Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Handle pandas timestamp with nanosecs precision #49370

Merged
merged 9 commits into from
Jan 4, 2025

Conversation

srinathk10
Copy link
Contributor

Why are these changes needed?

Handle pandas timestamp with nanosecs precision

Related issue number

"Closes #49297"

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@srinathk10 srinathk10 requested a review from a team as a code owner December 19, 2024 21:54
Copy link
Contributor

@omatthew98 omatthew98 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Few comments about the tests, I also wonder if we could somehow fold all these tests into a single parameterized test that is agnostic of the type (e.g. use the expected type in the parameters somewhere to know what type to expect. Would help with the standardization across the tests.

# Convert Python datetime to pandas Timestamp with nanosecond precision
value = pd.Timestamp(value)
value = pa.array([value], type=pa.timestamp("ns"))[0]

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are there any ways that this conversion can fail or precision can be lost? I imagine probably not but might be good to note anything here if there are.

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hey again, piping in from the original issue.

If this is already a datetime object, hasn't it already lost nanosecond precision? Is it too late to perform this coercion?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Default behavior is type coercion as we discovered in the bug from the test case. This is explicitly handling the type conversion for nanoseconds.

"df, expected_df",
[
(
create_timestamp_dataframe(),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Unless I am missing something this looks identical to the result of create_timestamp_dataframe? Should we just call that function once? Should we only pass in one dataframe?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah I see now that there is a 1ns difference, can we include a comment to specify that? Or could just have one DF passed in and do the addition of 1ns in the test itself?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I will check if I can simplify this without limiting the correctness check.

processed_df = result.to_pandas()
assert processed_df.shape == df.shape, "DataFrame shapes do not match"
pd.testing.assert_frame_equal(processed_df, expected_df)
assert (processed_df["timestamp"] - df["timestamp"]).iloc[0] == pd.Timedelta(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Are these last three asserts redundant with the above equality check?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ah this is checking for all 3 rows.

result = ray_data.map(process_timestamp_data)
processed_df = result.to_pandas()
# Ensure numpy.datetime64 is correctly converted to pandas Timestamp
assert isinstance(processed_df["timestamp"].iloc[0], pd.Timestamp)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we just check if the series for the column has the correct type?

assert isinstance(processed_df["timestamp"].iloc[0], pd.Timestamp)

# Check that the timestamp has been incremented by 1ns
assert (processed_df["timestamp"] - df["timestamp"]).min() == pd.Timedelta(1, "ns")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we standardize how we are checking the diff of 1ns? Seems like we are doing this, the equality check with the expected check and also individually checking the differences. Not super opinionated on which is the clearest but let's just use one (unless I am missing some subtlety here)
.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This one is datetime vs np.datetime64. Let me consolidate this code better.

@srinathk10 srinathk10 added the go add ONLY when ready to merge, run all tests label Dec 23, 2024
Copy link
Contributor

@omatthew98 omatthew98 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for cleaning up the tests! We could probably consolidate the two slightly more but approving regardless to unblock.

)
],
)
def test_map_numpy_datetime(df, expected_df, ray_start_regular_shared):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: It seems like test_map_numpy_datetime and test_map_timestamp_nanosecs have the same body? Can probably move this to be one test with two different sets of parameters.

if isinstance(value, (pd.Timestamp, np.datetime64)):
# If it's a pandas Timestamp or numpy datetime64, convert to pyarrow
# Timestamp
value = pa.array([value], type=pa.timestamp("ns"))[0]
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if the target block type is pandas, will converting it to pyarrow Timestamp be compatible?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should probably put this in the arrow_block.py subclass

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also, can you comment that the purpose of this conversion is to avoid losing precision?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if the target block type is pandas, will converting it to pyarrow Timestamp be compatible?

Here in add API, we are preventing implicit type coercion when we handing Pandas timestamp (nanoseconds) type while converting to Pyarrow Table.

Given Pyarrow is holding precision in Nanoseconds, converting Pyarrow -> Pandas does retain nanoseconds precision. Pytests convert the returned resultset (ds.take_all()) to Pandas and does verification of timestamp nanosecs as well.

)
],
)
def test_map_timestamp_nanosecs(df, expected_df, ray_start_regular_shared):
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • can you document the purpose of each test?
  • maybe put them in test_pandas.py, as the issue is specific to Pandas.

@srinathk10 srinathk10 force-pushed the srinathk10-timestamp-nanosecs branch from 229aeb1 to d63bf5f Compare January 3, 2025 02:51
Signed-off-by: Srinath Krishnamachari <[email protected]>
Signed-off-by: Srinath Krishnamachari <[email protected]>
Signed-off-by: Srinath Krishnamachari <[email protected]>
dtype=f"datetime64[{precision}]",
)
# Manually handle nanoseconds if the precision is 'ns'
def convert_to_datetime64(dt: datetime) -> np.datetime64:
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit, would be cleaner to define it as a standalone function

pandas_builder.add(row)
pandas_block = pandas_builder.build()

# assert pd.api.types.is_datetime64_ns_dtype(pandas_block["col2"])
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is it still needed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oops this needs to be uncommented.

@@ -45,28 +45,76 @@ def validate_numpy_batch(batch: Union[Dict[str, np.ndarray], Dict[str, list]]) -


def _detect_highest_datetime_precision(datetime_list: List[datetime]) -> str:
"""Detect the highest precision for a list of datetime objects.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

didn't know there is such a function. This looks nicer than the previous approach. 👍

Copy link
Contributor

@omatthew98 omatthew98 left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Two comments to clean up the _convert_datetime_list_to_array function.

# Manually calculate nanoseconds by adding microseconds * 1000
nanoseconds = dt.microsecond * 1000 + dt.nanosecond
# Now manually create a datetime64 with nanosecond precision
return np.datetime64(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I wonder if for this case, we could simply add a timedelta to the lower precision np.datetime created from the original time stamp. This would prevent us from having to format the string directly, which is probably fine. I am envisioning something like this (from ChatGPT):

import datetime
import numpy as np

# Suppose we have year=2025, month=1, day=2, hour=15, minute=30, second=45,
# plus an extra 789 nanoseconds beyond microseconds.
# 1 microsecond = 1000 nanoseconds
# So total ns = (123456 * 1000) + 789
# But standard datetime can only store the microseconds part (123456):
base_dt = datetime.datetime(2025, 1, 2, 15, 30, 45, 123456)
leftover_ns = 789

# Convert the base datetime to a ns-based np.datetime64
base_np_dt = np.datetime64(base_dt, 'ns')

# Add the leftover nanoseconds as a timedelta64:
final_np_dt = base_np_dt + np.timedelta64(leftover_ns, 'ns')

print(final_np_dt)
# 2025-01-02T15:30:45.123456789

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nice will give this a shot. Yea, converting to string was not ideal, but I was trying a lot of options and none worked so far.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah from what I read it seems like unless the above works (I didn't do too much manual testing) the string formatting is unfortunately what we have to go with. Seems fine to leave as I don't think we are going to be editing this code much, but could be nice to go with the above.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This worked :).

f"{dt.second:02d}.{nanoseconds:09d}",
"ns",
)
elif precision == "us":
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Nit: Can collapse a lot / all of these into the same branch with something like return np.datetime64(dt).astype(f"datetime64[{precision}]")

Signed-off-by: Srinath Krishnamachari <[email protected]>
Signed-off-by: Srinath Krishnamachari <[email protected]>
Signed-off-by: Srinath Krishnamachari <[email protected]>
Signed-off-by: Srinath Krishnamachari <[email protected]>
@raulchen raulchen merged commit 3df8163 into master Jan 4, 2025
5 checks passed
@raulchen raulchen deleted the srinathk10-timestamp-nanosecs branch January 4, 2025 18:21
roshankathawate pushed a commit to roshankathawate/ray that referenced this pull request Jan 7, 2025
## Why are these changes needed?
Handle pandas timestamp with nanosecs precision

## Related issue number

"Closes ray-project#49297"

---------

Signed-off-by: Srinath Krishnamachari <[email protected]>
roshankathawate pushed a commit to roshankathawate/ray that referenced this pull request Jan 9, 2025
## Why are these changes needed?
Handle pandas timestamp with nanosecs precision

## Related issue number

"Closes ray-project#49297"

---------

Signed-off-by: Srinath Krishnamachari <[email protected]>
Signed-off-by: Roshan Kathawate <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

[Data] Inconsistent behavior with Ray Data and timestamps
4 participants